Skip to content

KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module#13049

Merged
ijuma merged 7 commits intoapache:trunkfrom
ijuma:kafka-14478-move-log-config-storage
Jan 4, 2023
Merged

KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module#13049
ijuma merged 7 commits intoapache:trunkfrom
ijuma:kafka-14478-move-log-config-storage

Conversation

@ijuma
Copy link
Member

@ijuma ijuma commented Dec 28, 2022

Additional notable changes to fix multiple dependency ordering issues:

  • Moved ConfigSynonym to server-common
  • Moved synonyms from LogConfig to ServerTopicConfigSynonyms
  • Removed LogConfigDef define overrides and rely on
    ServerTopicConfigSynonyms instead.
  • Moved LogConfig.extractLogConfigMap to KafkaConfig
  • Consolidated relevant defaults from KafkaConfig/LogConfig in the latter
  • Consolidate relevant config name definitions in TopicConfig
  • Move ThrottledReplicaListValidator to storage

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@ijuma ijuma changed the title KAFKA-14478: Move LogConfig and related to storage module KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module Dec 28, 2022
@ijuma ijuma force-pushed the kafka-14478-move-log-config-storage branch from 9dbdb87 to 9532d07 Compare December 28, 2022 02:55
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed the trim from here since we do it in ensureValid as well.

@ijuma ijuma force-pushed the kafka-14478-move-log-config-storage branch 3 times, most recently from aaed51e to 4277ed7 Compare December 28, 2022 14:56
ijuma added a commit that referenced this pull request Dec 28, 2022
…MetadataLog` (#13051)

`LogConfig.validateValues` may fail or incorrectly succeed if the properties don't include defaults.

During the conversion of `LogConfig` to Java (#13049), it became clear that the `asInstanceOf[Long]`
calls in `LogConfig.validateValues` were converting `null` to `0` when this method was invoked
from `KafkaMetadataLog`. This means that it would be possible for it to validate successfully
in cases where it should not.

Reviewers: José Armando García Sancio <jsancio@apache.org>
@ijuma ijuma force-pushed the kafka-14478-move-log-config-storage branch from 6bf3b84 to f5946bd Compare December 28, 2022 18:56
maxBatchSizeInBytes: Int,
maxFetchSizeInBytes: Int,
fileDeleteDelayMs: Int,
fileDeleteDelayMs: Long,
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed to long to match the type of the config (even if the default value happened to fit an int before).

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch!

.define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc)
.define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc)
.define(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc)
.define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc)
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's a bit unintuitive that LogFlushSchedulerIntervalMsProp (vs LogFlushIntervalMsProp) defaults to LogConfig.DEFAULT_FLUSH_MS, but this preserves the existing approach. There is logic to set LogFlushIntervalMsProp to LogFlushSchedulerIntervalMsProp if unset.

@ijuma
Copy link
Member Author

ijuma commented Dec 28, 2022

JDK 8 and JDK 11 builds passed, JDK 17 had one unrelated failure.

if (segmentJitterMs == 0)
return 0;
else
return Utils.abs(ThreadLocalRandom.current().nextInt()) % Math.min(segmentJitterMs, segmentMs);
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Slight change here: we now use a ThreadLocalRandom instead of a global Random.

@ijuma ijuma marked this pull request as ready for review December 28, 2022 21:48
@ijuma ijuma requested review from junrao and satishd December 28, 2022 21:55
TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG,
TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG,
LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG,
FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG and FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG to this list since they never had server defaults.

}

// Visible for testing
public static class LogConfigDef extends ConfigDef {
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed define overrides since we now rely on the statically defined ServerTopicConfigSynonyms to retrieve the server config default/synonym.

@ijuma
Copy link
Member Author

ijuma commented Jan 2, 2023

@satishd If you have cycles, this PR is ready for review.

Copy link
Member

@mimaison mimaison left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. I left a few small suggestions.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe put it with the other java.util imports below

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could inline these now

"all replicas for this topic.";

@SuppressWarnings("deprecation")
private static final String MESSAGE_FORMAT_VERSION_DOC = TopicConfig.MESSAGE_FORMAT_VERSION_DOC;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need this constant? We already suppress deprecation in the static block below.

Copy link
Member Author

@ijuma ijuma Jan 3, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do, there's no way to suppress deprecations within the static block (I made this change to get the code to compile). If you find a way to avoid this constant, please do share.

private final int maxMessageSize;
private final Map<?, ?> props;

public LogConfig(Map<?, ?> props) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Considering the number of callers with new Properties() in tests I wonder if we should have a zero argument constructor too.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would prefer to leave it as is. Generally, I don't think it's an improvement overall to add overloads that are only used in tests.

@ijuma
Copy link
Member Author

ijuma commented Jan 3, 2023

@mimaison Thanks for the review. I addressed or replied to each comment.

Copy link
Member

@satishd satishd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @ijuma for the PR. LGTM

@ijuma ijuma merged commit 96d9710 into apache:trunk Jan 4, 2023
@ijuma ijuma deleted the kafka-14478-move-log-config-storage branch January 4, 2023 10:42
@ijuma
Copy link
Member Author

ijuma commented Jan 4, 2023

I went ahead and merged this to unblock other PRs, but happy to follow up if @mimaison or @junrao have additional comments.

Copy link
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ijuma : Thanks for the PR. Just a couple of minor comments.

validateValues(valueMaps);
}

public static void main(String[] args) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we moved the package of this tool, should we change the following line in build.gradle?

mainClass = 'kafka.log.LogConfig'

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I observed this issue when I pulled the trunk changes in other PR and raised a fix for that in #13079

return Utils.mkEntry(topicConfigName, synonymsWithPrefix);
}

private static Entry<String, List<ConfigSynonym>> single(String topicConfigName, String brokerConfigName) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would differentName be more intuitive than single?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good question. The naming scheme was trying to cover 3 different cases, each with prefix/no prefix variants:

  1. Single mapping between topic and broker config name: single (this is like Collections.singleton in a way)
  2. Mapping of topic to multiple broker synonyms: list
  3. A mapping where the topic and broker config have the same name: sameName (this could also have been called identity now that I think of it).

Are you suggesting that all single* methods become differentName*? What about list*, does that become differentNameList*?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the explanation. Got it. I am not sure that I could make the naming better. We can just keep it as it is.

guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…MetadataLog` (apache#13051)

`LogConfig.validateValues` may fail or incorrectly succeed if the properties don't include defaults.

During the conversion of `LogConfig` to Java (apache#13049), it became clear that the `asInstanceOf[Long]`
calls in `LogConfig.validateValues` were converting `null` to `0` when this method was invoked
from `KafkaMetadataLog`. This means that it would be possible for it to validate successfully
in cases where it should not.

Reviewers: José Armando García Sancio <jsancio@apache.org>
guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…le (apache#13049)

Additional notable changes to fix multiple dependency ordering issues:

* Moved `ConfigSynonym` to `server-common`
* Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms `
* Removed `LogConfigDef` `define` overrides and rely on
   `ServerTopicConfigSynonyms` instead.
* Moved `LogConfig.extractLogConfigMap` to `KafkaConfig`
* Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter
* Consolidate relevant config name definitions in `TopicConfig`
* Move `ThrottledReplicaListValidator` to `storage`

Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants

Comments